79868b
@@ -10,6 +10,7 @@
  */
 package org.apache.hadoop.hbase.replication.master;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
@@ -41,12 +42,11 @@
import org.apache.zookeeper.KeeperException;
  * deleting it from hfile archive directory.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate implements Abortable {
+public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
   private static final Log LOG = LogFactory.getLog(ReplicationHFileCleaner.class);
   private ZooKeeperWatcher zkw;
   private ReplicationQueuesClient rqc;
   private boolean stopped = false;
-  private boolean aborted;
 
   @Override
   public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
@@ -129,18 +129,27 @@
public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate implements
     // Make my own Configuration. Then I'll have my own connection to zk that
     // I can close myself when time comes.
     Configuration conf = new Configuration(config);
+    try {
+      setConf(conf, new ZooKeeperWatcher(conf, "replicationHFileCleaner", null));
+    } catch (IOException e) {
+      LOG.error("Error while configuring " + this.getClass().getName(), e);
+    }
+  }
+
+  @VisibleForTesting
+  public void setConf(Configuration conf, ZooKeeperWatcher zk) {
     super.setConf(conf);
     try {
-      initReplicationQueuesClient(conf);
+      initReplicationQueuesClient(conf, zk);
     } catch (IOException e) {
       LOG.error("Error while configuring " + this.getClass().getName(), e);
     }
   }
 
-  private void initReplicationQueuesClient(Configuration conf)
+  private void initReplicationQueuesClient(Configuration conf, ZooKeeperWatcher zk)
       throws ZooKeeperConnectionException, IOException {
-    this.zkw = new ZooKeeperWatcher(conf, "replicationHFileCleaner", null);
-    this.rqc = ReplicationFactory.getReplicationQueuesClient(zkw, conf, this);
+    this.zkw = zk;
+    this.rqc = ReplicationFactory.getReplicationQueuesClient(zkw, conf, new WarnOnlyAbortable());
   }
 
   @Override
@@ -160,18 +169,6 @@
public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate implements
     return this.stopped;
   }
 
-  @Override
-  public void abort(String why, Throwable e) {
-    LOG.warn("Aborting ReplicationHFileCleaner because " + why, e);
-    this.aborted = true;
-    stop(why);
-  }
-
-  @Override
-  public boolean isAborted() {
-    return this.aborted;
-  }
-
   @Override
   public boolean isFileDeletable(FileStatus fStat) {
     Set<String> hfileRefsFromQueue;
@@ -190,4 +187,19 @@
public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate implements
     }
     return !hfileRefsFromQueue.contains(fStat.getPath().getName());
   }
+
+  private static class WarnOnlyAbortable implements Abortable {
+    @Override
+    public void abort(String why, Throwable e) {
+      LOG.warn("ReplicationHFileCleaner received abort, ignoring.  Reason: " + why);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(e);
+      }
+    }
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+  }
 }
